diff --git a/.build/cassandra-deps-template.xml b/.build/cassandra-deps-template.xml index b7b38afddb34..47e8a6cda676 100644 --- a/.build/cassandra-deps-template.xml +++ b/.build/cassandra-deps-template.xml @@ -376,5 +376,9 @@ io.github.jbellis jvector + + com.vdurmont + semver4j + diff --git a/.build/parent-pom-template.xml b/.build/parent-pom-template.xml index 3991bc5fdeac..54e373f42afc 100644 --- a/.build/parent-pom-template.xml +++ b/.build/parent-pom-template.xml @@ -1227,6 +1227,11 @@ big-math 2.3.0 + + com.vdurmont + semver4j + 3.1.0 + diff --git a/CHANGES.txt b/CHANGES.txt index 940523336503..d90ee5de251b 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,4 +1,5 @@ 5.0-beta2 + * Add a startup check to fail startup when using invalid configuration with certain Kernel and FS type (CASSANDRA-19196) * UCS min_sstable_size should not be lower than target_sstable_size lower bound (CASSANDRA-19112) * Fix the correspondingMessagingVersion of SSTable format and improve TTL overflow tests coverage (CASSANDRA-19197) * Fix resource cleanup after SAI query timeouts (CASSANDRA-19177) diff --git a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java index 53341c4c5fba..16a24fd2f01c 100644 --- a/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java +++ b/src/java/org/apache/cassandra/config/CassandraRelevantProperties.java @@ -264,6 +264,9 @@ public enum CassandraRelevantProperties @Deprecated(since = "4.1") IGNORE_DC("cassandra.ignore_dc"), IGNORE_DYNAMIC_SNITCH_SEVERITY("cassandra.ignore_dynamic_snitch_severity"), + + IGNORE_KERNEL_BUG_1057843_CHECK("cassandra.ignore_kernel_bug_1057843_check"), + IGNORE_MISSING_NATIVE_FILE_HINTS("cassandra.require_native_file_hints"), /** @deprecated should be removed in favor of enable flag of relevant startup check (checkRack) */ /** @deprecated See CASSANDRA-17797 */ diff --git a/src/java/org/apache/cassandra/config/Config.java b/src/java/org/apache/cassandra/config/Config.java index a4dcae506448..0299fa348451 100644 --- a/src/java/org/apache/cassandra/config/Config.java +++ b/src/java/org/apache/cassandra/config/Config.java @@ -1147,7 +1147,12 @@ public enum DiskAccessMode mmap_index_only, standard, legacy, - direct // Direct-I/O is enabled for commitlog disk only. + + /** + * Direct-I/O is enabled for commitlog disk only. + * When adding support for direct IO, update {@link org.apache.cassandra.service.StartupChecks#checkKernelBug1057843} + */ + direct } public enum MemtableAllocationType diff --git a/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java b/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java index d27a93555b76..3e157b6b9a01 100644 --- a/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java +++ b/src/java/org/apache/cassandra/io/util/DataOutputBuffer.java @@ -21,11 +21,14 @@ import java.nio.BufferOverflowException; import java.nio.ByteBuffer; import java.nio.channels.WritableByteChannel; +import java.nio.charset.CharacterCodingException; +import java.nio.charset.StandardCharsets; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import io.netty.util.concurrent.FastThreadLocal; +import org.apache.cassandra.utils.ByteBufferUtil; import static org.apache.cassandra.config.CassandraRelevantProperties.DATA_OUTPUT_BUFFER_ALLOCATE_TYPE; import static org.apache.cassandra.config.CassandraRelevantProperties.DOB_DOUBLING_THRESHOLD_MB; @@ -286,4 +289,16 @@ public byte[] toByteArray() buffer.get(result); return result; } + + public String asString() + { + try + { + return ByteBufferUtil.string(buffer(), StandardCharsets.UTF_8); + } + catch (CharacterCodingException e) + { + throw new RuntimeException(e); + } + } } diff --git a/src/java/org/apache/cassandra/service/StartupChecks.java b/src/java/org/apache/cassandra/service/StartupChecks.java index d04d559a3bcd..934a17b06494 100644 --- a/src/java/org/apache/cassandra/service/StartupChecks.java +++ b/src/java/org/apache/cassandra/service/StartupChecks.java @@ -44,10 +44,12 @@ import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; +import com.google.common.collect.Range; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.vdurmont.semver4j.Semver; import net.jpountz.lz4.LZ4Factory; import org.apache.cassandra.config.CassandraRelevantProperties; import org.apache.cassandra.config.Config; @@ -75,6 +77,7 @@ import static org.apache.cassandra.config.CassandraRelevantProperties.CASSANDRA_JMX_LOCAL_PORT; import static org.apache.cassandra.config.CassandraRelevantProperties.COM_SUN_MANAGEMENT_JMXREMOTE_PORT; +import static org.apache.cassandra.config.CassandraRelevantProperties.IGNORE_KERNEL_BUG_1057843_CHECK; import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VERSION; import static org.apache.cassandra.config.CassandraRelevantProperties.JAVA_VM_NAME; import static org.apache.cassandra.utils.Clock.Global.currentTimeMillis; @@ -129,7 +132,8 @@ public enum StartupCheckType // The default set of pre-flight checks to run. Order is somewhat significant in that we probably // always want the system keyspace check run last, as this actually loads the schema for that // keyspace. All other checks should not require any schema initialization. - private final List DEFAULT_TESTS = ImmutableList.of(checkJemalloc, + private final List DEFAULT_TESTS = ImmutableList.of(checkKernelBug1057843, + checkJemalloc, checkLz4Native, checkValidLaunchDate, checkJMXPorts, @@ -187,6 +191,64 @@ public void verify(StartupChecksOptions options) throws StartupException } } + // https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=1057843 + public static final StartupCheck checkKernelBug1057843 = new StartupCheck() + { + @Override + public void execute(StartupChecksOptions startupChecksOptions) throws StartupException + { + if (startupChecksOptions.isDisabled(getStartupCheckType())) + return; + + if (!FBUtilities.isLinux) + return; + + Set directIOWritePaths = new HashSet<>(); + if (DatabaseDescriptor.getCommitLogWriteDiskAccessMode() == Config.DiskAccessMode.direct) + directIOWritePaths.add(new File(DatabaseDescriptor.getCommitLogLocation()).toPath()); + // TODO: add data directories when direct IO is supported for flushing and compaction + + if (!directIOWritePaths.isEmpty() && IGNORE_KERNEL_BUG_1057843_CHECK.getBoolean()) + { + logger.info("Ignoring check for the kernel bug 1057843 against the following paths configured to be accessed with Direct IO: {}", directIOWritePaths); + return; + } + + Set affectedFileSystemTypes = Set.of("ext4"); + Set affectedPaths = new HashSet<>(); + for (Path path : directIOWritePaths) + { + try + { + if (affectedFileSystemTypes.contains(Files.getFileStore(path).type().toLowerCase())) + affectedPaths.add(path); + } + catch (IOException e) + { + throw new StartupException(StartupException.ERR_WRONG_MACHINE_STATE, "Failed to determine file system type for path " + path, e); + } + } + + if (affectedPaths.isEmpty()) + return; + + Range affectedKernels = Range.closedOpen(new Semver("6.1.64", Semver.SemverType.LOOSE), + new Semver("6.1.66", Semver.SemverType.LOOSE)); + + Semver kernelVersion = FBUtilities.getKernelVersion(); + if (!affectedKernels.contains(kernelVersion.withClearedSuffixAndBuild())) + return; + + throw new StartupException(StartupException.ERR_WRONG_MACHINE_STATE, + String.format("Detected kernel version %s with affected file system types %s and direct IO enabled for paths %s. " + + "This combination is known to cause data corruption. To start Cassandra in this environment, " + + "you have to disable direct IO for the affected paths. If you are sure the verification provided " + + "a false positive result, you can suppress it by setting '" + IGNORE_KERNEL_BUG_1057843_CHECK.getKey() + "' system property to 'true'. " + + "Please see https://bugs.debian.org/cgi-bin/bugreport.cgi?bug=1057843 for more information.", + kernelVersion, affectedFileSystemTypes, affectedPaths)); + } + }; + public static final StartupCheck checkJemalloc = new StartupCheck() { @Override diff --git a/src/java/org/apache/cassandra/utils/FBUtilities.java b/src/java/org/apache/cassandra/utils/FBUtilities.java index fe31ecd3a628..444864ddc7b4 100644 --- a/src/java/org/apache/cassandra/utils/FBUtilities.java +++ b/src/java/org/apache/cassandra/utils/FBUtilities.java @@ -34,8 +34,10 @@ import java.nio.ByteBuffer; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; +import java.time.Duration; import java.time.Instant; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.Comparator; @@ -45,11 +47,13 @@ import java.util.NavigableSet; import java.util.Optional; import java.util.Properties; +import java.util.Scanner; import java.util.TreeSet; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.function.Supplier; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.zip.CRC32; @@ -59,11 +63,14 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; +import com.google.common.base.Suppliers; import com.google.common.collect.ImmutableList; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.vdurmont.semver4j.Semver; import org.apache.cassandra.audit.IAuditLogger; import org.apache.cassandra.auth.AllowAllNetworkAuthorizer; import org.apache.cassandra.auth.IAuthenticator; @@ -131,11 +138,19 @@ public class FBUtilities private static int availableProcessors = CASSANDRA_AVAILABLE_PROCESSORS.getInt(DatabaseDescriptor.getAvailableProcessors()); + private static volatile Supplier kernelVersionSupplier = Suppliers.memoize(FBUtilities::getKernelVersionFromUname); + public static void setAvailableProcessors(int value) { availableProcessors = value; } + @VisibleForTesting + public static void setKernelVersionSupplier(Supplier supplier) + { + kernelVersionSupplier = supplier; + } + public static int getAvailableProcessors() { if (availableProcessors > 0) @@ -1080,6 +1095,66 @@ public static void exec(ProcessBuilder pb) throws IOException } } + /** + * Starts and waits for the given cmd to finish. If the process does not finish within timeout, + * it will be destroyed. + * + * @param env additional environment variables to set + * @param timeout timeout for the process to finish, or zero/null to wait forever + * @param outBufSize the maximum size of the collected std output; the overflow will be discarded + * @param errBufSize the maximum size of the collected std error; the overflow will be discarded + * @param cmd the command to execute + * @return the std output of the process up to the size specified by outBufSize + */ + public static String exec(Map env, Duration timeout, int outBufSize, int errBufSize, String... cmd) throws IOException, TimeoutException, InterruptedException + { + if (env == null) + env = Map.of(); + if (timeout == null) + timeout = Duration.ZERO; + + ProcessBuilder processBuilder = new ProcessBuilder(cmd); + processBuilder.environment().putAll(env); + Process process = processBuilder.start(); + try (DataOutputBuffer err = new DataOutputBuffer(); + DataOutputBuffer out = new DataOutputBuffer(); + OutputStream overflowSink = OutputStream.nullOutputStream()) + { + boolean completed; + if (timeout.isZero()) + { + process.waitFor(); + completed = true; + } + else + { + completed = process.waitFor(timeout.toMillis(), TimeUnit.MILLISECONDS); + } + + copy(process.getInputStream(), out, outBufSize); + long outOverflow = process.getInputStream().transferTo(overflowSink); + + copy(process.getErrorStream(), err, errBufSize); + long errOverflow = process.getErrorStream().transferTo(overflowSink); + + if (!completed) + { + process.destroyForcibly(); + logger.error("Command {} did not complete in {}, killed forcibly:\noutput:\n{}\n(truncated {} bytes)\nerror:\n{}\n(truncated {} bytes)", + Arrays.toString(cmd), timeout, out.asString(), outOverflow, err.asString(), errOverflow); + throw new TimeoutException("Command " + Arrays.toString(cmd) + " did not complete in " + timeout); + } + int r = process.exitValue(); + if (r != 0) + { + logger.error("Command {} failed with exit code {}:\noutput:\n{}\n(truncated {} bytes)\nerror:\n{}\n(truncated {} bytes)", + Arrays.toString(cmd), r, out.asString(), outOverflow, err.asString(), errOverflow); + throw new IOException("Command " + Arrays.toString(cmd) + " failed with exit code " + r); + } + return out.asString(); + } + } + public static void updateChecksumInt(Checksum checksum, int v) { checksum.update((v >>> 24) & 0xFF); @@ -1320,4 +1395,53 @@ public static void closeQuietly(Object o) logger.warn("Closing {} had an unexpected exception", o, e); } } + + public static Semver getKernelVersion() + { + return kernelVersionSupplier.get(); + } + + @VisibleForTesting + static Semver getKernelVersionFromUname() + { + // TODO rewrite this method with Oshi when it is eventually included in the project + if (!isLinux) + return null; + + try + { + String output = exec(Map.of(), Duration.ofSeconds(5), 1024, 1024, "uname", "-r"); + + if (output.isEmpty()) + throw new RuntimeException("Error while trying to get kernel version, 'uname -r' returned empty output"); + + return parseKernelVersion(output); + } + catch (IOException | TimeoutException e) + { + throw new RuntimeException("Error while trying to get kernel version", e); + } + catch (InterruptedException e) + { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + + @VisibleForTesting + static Semver parseKernelVersion(String versionString) + { + Preconditions.checkNotNull(versionString, "kernel version cannot be null"); + try (Scanner scanner = new Scanner(versionString)) + { + while (scanner.hasNextLine()) + { + String version = scanner.nextLine().trim(); + if (version.isEmpty()) + continue; + return new Semver(version, Semver.SemverType.LOOSE); + } + } + throw new IllegalArgumentException("Error while trying to parse kernel version - no version found"); + } } \ No newline at end of file diff --git a/test/unit/org/apache/cassandra/service/StartupChecksTest.java b/test/unit/org/apache/cassandra/service/StartupChecksTest.java index 6a26899beb8f..6fb063987ad0 100644 --- a/test/unit/org/apache/cassandra/service/StartupChecksTest.java +++ b/test/unit/org/apache/cassandra/service/StartupChecksTest.java @@ -18,32 +18,57 @@ package org.apache.cassandra.service; import java.io.IOException; +import java.nio.file.FileStore; +import java.nio.file.FileSystem; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.nio.file.spi.FileSystemProvider; import java.time.Instant; import java.util.List; - -import org.apache.cassandra.config.StartupChecksOptions; -import org.apache.cassandra.io.util.File; -import org.junit.*; - +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.stream.Collectors; + +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +import com.vdurmont.semver4j.Semver; import org.apache.cassandra.SchemaLoader; +import org.apache.cassandra.config.Config.DiskAccessMode; import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.schema.SchemaConstants; -import org.apache.cassandra.db.*; +import org.apache.cassandra.config.StartupChecksOptions; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Directories; +import org.apache.cassandra.db.Keyspace; import org.apache.cassandra.exceptions.StartupException; +import org.apache.cassandra.io.filesystem.ForwardingFileSystem; +import org.apache.cassandra.io.filesystem.ForwardingFileSystemProvider; +import org.apache.cassandra.io.filesystem.ForwardingPath; +import org.apache.cassandra.io.util.File; +import org.apache.cassandra.schema.SchemaConstants; import org.apache.cassandra.service.DataResurrectionCheck.Heartbeat; import org.apache.cassandra.utils.Clock; +import org.apache.cassandra.utils.FBUtilities; import static java.util.Collections.singletonList; import static org.apache.cassandra.config.CassandraRelevantProperties.TEST_INVALID_LEGACY_SSTABLE_ROOT; import static org.apache.cassandra.io.util.FileUtils.createTempFile; import static org.apache.cassandra.service.DataResurrectionCheck.HEARTBEAT_FILE_CONFIG_PROPERTY; import static org.apache.cassandra.service.StartupChecks.StartupCheckType.check_data_resurrection; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class StartupChecksTest { @@ -204,6 +229,104 @@ List getTablesGcPeriods(String userKeyspace) verifyFailure(startupChecks, "Invalid tables: abc.def"); } + @Test + public void testKernelBug1057843Check() throws Exception + { + Assume.assumeTrue(DatabaseDescriptor.getCommitLogCompression() == null); // we would not be able to enable direct io otherwise + testKernelBug1057843Check("ext4", DiskAccessMode.direct, new Semver("6.1.63.1-generic"), false); + testKernelBug1057843Check("ext4", DiskAccessMode.direct, new Semver("6.1.64.1-generic"), true); + testKernelBug1057843Check("ext4", DiskAccessMode.direct, new Semver("6.1.65.1-generic"), true); + testKernelBug1057843Check("ext4", DiskAccessMode.direct, new Semver("6.1.66.1-generic"), false); + testKernelBug1057843Check("tmpfs", DiskAccessMode.direct, new Semver("6.1.64.1-generic"), false); + testKernelBug1057843Check("ext4", DiskAccessMode.mmap, new Semver("6.1.64.1-generic"), false); + } + + private void withPathOverriddingFileSystem(Map pathOverrides, Callable callable) throws Exception + { + Map fileStores = Set.copyOf(pathOverrides.values()).stream().collect(Collectors.toMap(s -> s, s -> { + FileStore fs = mock(FileStore.class); + when(fs.type()).thenReturn(s); + return fs; + })); + FileSystem savedFileSystem = File.unsafeGetFilesystem(); + try + { + ForwardingFileSystemProvider fsp = new ForwardingFileSystemProvider(savedFileSystem.provider()) + { + @Override + public FileStore getFileStore(Path path) throws IOException + { + String override = pathOverrides.get(path.toString()); + if (override != null) + return fileStores.get(override); + + return super.getFileStore(path); + } + }; + + ForwardingFileSystem fs = new ForwardingFileSystem(File.unsafeGetFilesystem()) + { + private final FileSystem thisFileSystem = this; + + @Override + public FileSystemProvider provider() + { + return fsp; + } + + @Override + protected Path wrap(Path p) + { + return new ForwardingPath(p) + { + @Override + public FileSystem getFileSystem() + { + return thisFileSystem; + } + }; + } + }; + File.unsafeSetFilesystem(fs); + callable.call(); + } + finally + { + File.unsafeSetFilesystem(savedFileSystem); + } + } + + private void testKernelBug1057843Check(String fsType, DiskAccessMode diskAccessMode, Semver kernelVersion, boolean expectToFail) throws Exception + { + String commitLogLocation = Files.createTempDirectory("testKernelBugCheck").toString(); + + String savedCommitLogLocation = DatabaseDescriptor.getCommitLogLocation(); + DiskAccessMode savedCommitLogWriteDiskAccessMode = DatabaseDescriptor.getCommitLogWriteDiskAccessMode(); + Semver savedKernelVersion = FBUtilities.getKernelVersion(); + try + { + DatabaseDescriptor.setCommitLogLocation(commitLogLocation); + DatabaseDescriptor.setCommitLogWriteDiskAccessMode(diskAccessMode); + DatabaseDescriptor.initializeCommitLogDiskAccessMode(); + assertThat(DatabaseDescriptor.getCommitLogWriteDiskAccessMode()).isEqualTo(diskAccessMode); + FBUtilities.setKernelVersionSupplier(() -> kernelVersion); + withPathOverriddingFileSystem(Map.of(commitLogLocation, fsType), () -> { + if (expectToFail) + assertThatExceptionOfType(StartupException.class).isThrownBy(() -> StartupChecks.checkKernelBug1057843.execute(options)); + else + StartupChecks.checkKernelBug1057843.execute(options); + return null; + }); + } + finally + { + DatabaseDescriptor.setCommitLogLocation(savedCommitLogLocation); + DatabaseDescriptor.setCommitLogWriteDiskAccessMode(savedCommitLogWriteDiskAccessMode); + DatabaseDescriptor.initializeCommitLogDiskAccessMode(); + FBUtilities.setKernelVersionSupplier(() -> savedKernelVersion); + } + } + private void copyInvalidLegacySSTables(Path targetDir) throws IOException { File legacySSTableRoot = new File(Paths.get(TEST_INVALID_LEGACY_SSTABLE_ROOT.getString(), diff --git a/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java b/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java index cf54a6223b14..7b2bd88afdb8 100644 --- a/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java +++ b/test/unit/org/apache/cassandra/utils/FBUtilitiesTest.java @@ -23,13 +23,13 @@ import java.nio.ByteBuffer; import java.nio.charset.CharacterCodingException; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Arrays; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Random; import java.util.TreeMap; -import java.util.ArrayList; -import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -38,20 +38,30 @@ import java.util.concurrent.TimeUnit; import com.google.common.primitives.Ints; - import org.junit.Assert; +import org.junit.Assume; import org.junit.Test; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.cassandra.db.marshal.*; -import org.apache.cassandra.dht.*; - +import com.vdurmont.semver4j.Semver; import org.apache.cassandra.config.Config; import org.apache.cassandra.config.DatabaseDescriptor; - +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.ListType; +import org.apache.cassandra.db.marshal.UUIDType; +import org.apache.cassandra.dht.ByteOrderedPartitioner; +import org.apache.cassandra.dht.IPartitioner; +import org.apache.cassandra.dht.LengthPartitioner; +import org.apache.cassandra.dht.LocalPartitioner; +import org.apache.cassandra.dht.Murmur3Partitioner; +import org.apache.cassandra.dht.OrderPreservingPartitioner; +import org.apache.cassandra.dht.RandomPartitioner; + +import static org.apache.cassandra.utils.FBUtilities.parseKernelVersion; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -363,4 +373,27 @@ public void testPrettyPrintAndParseRandom() Assert.assertEquals(value, FBUtilities.parseHumanReadable(vDec, sep, unit), getDelta(value)); } } + + @Test + public void testParseKernelVersion() + { + assertThat(parseKernelVersion("4.4.0-21-generic").toString()).isEqualTo("4.4.0-21-generic"); + assertThat(parseKernelVersion("4.4.0-pre21-generic").toString()).isEqualTo("4.4.0-pre21-generic"); + assertThat(parseKernelVersion("4.4-pre21-generic").toString()).isEqualTo("4.4-pre21-generic"); + assertThat(parseKernelVersion("4.4.0-21-generic\n").toString()).isEqualTo("4.4.0-21-generic"); + assertThat(parseKernelVersion("\n4.4.0-21-generic\n").toString()).isEqualTo("4.4.0-21-generic"); + assertThat(parseKernelVersion("\n 4.4.0-21-generic \n").toString()).isEqualTo("4.4.0-21-generic"); + + assertThatExceptionOfType(IllegalArgumentException.class).isThrownBy(() -> parseKernelVersion("\n \n")) + .withMessageContaining("no version found"); + } + + @Test + public void testGetKernelVersion() + { + Assume.assumeTrue(FBUtilities.isLinux); + Semver kernelVersion = FBUtilities.getKernelVersion(); + assertThat(kernelVersion).isGreaterThan(new Semver("0.0.0", Semver.SemverType.LOOSE)); + assertThat(kernelVersion).isLessThan(new Semver("100.0.0", Semver.SemverType.LOOSE)); + } }