Skip to content

Commit

Permalink
Add a startup check to fail startup when using invalid configuration …
Browse files Browse the repository at this point in the history
…with certain Kernel and FS type

Patch by Jacek Lewandowski; reviewed by Maxwell Guo, Stefan Miklosovic for CASSANDRA-19196
  • Loading branch information
jacek-lewandowski committed Dec 21, 2023
1 parent 3b05051 commit f0ea12c
Show file tree
Hide file tree
Showing 10 changed files with 392 additions and 17 deletions.
4 changes: 4 additions & 0 deletions .build/cassandra-deps-template.xml
Expand Up @@ -376,5 +376,9 @@
<groupId>io.github.jbellis</groupId>
<artifactId>jvector</artifactId>
</dependency>
<dependency>
<groupId>com.vdurmont</groupId>
<artifactId>semver4j</artifactId>
</dependency>
</dependencies>
</project>
5 changes: 5 additions & 0 deletions .build/parent-pom-template.xml
Expand Up @@ -1227,6 +1227,11 @@
<artifactId>big-math</artifactId>
<version>2.3.0</version>
</dependency>
<dependency>
<groupId>com.vdurmont</groupId>
<artifactId>semver4j</artifactId>
<version>3.1.0</version>
</dependency>
</dependencies>
</dependencyManagement>
</project>
1 change: 1 addition & 0 deletions 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)
Expand Down
Expand Up @@ -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 */
Expand Down
7 changes: 6 additions & 1 deletion src/java/org/apache/cassandra/config/Config.java
Expand Up @@ -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
Expand Down
15 changes: 15 additions & 0 deletions src/java/org/apache/cassandra/io/util/DataOutputBuffer.java
Expand Up @@ -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;
Expand Down Expand Up @@ -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);
}
}
}
64 changes: 63 additions & 1 deletion src/java/org/apache/cassandra/service/StartupChecks.java
Expand Up @@ -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;
Expand Down Expand Up @@ -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;
Expand Down Expand Up @@ -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<StartupCheck> DEFAULT_TESTS = ImmutableList.of(checkJemalloc,
private final List<StartupCheck> DEFAULT_TESTS = ImmutableList.of(checkKernelBug1057843,
checkJemalloc,
checkLz4Native,
checkValidLaunchDate,
checkJMXPorts,
Expand Down Expand Up @@ -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<Path> 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<String> affectedFileSystemTypes = Set.of("ext4");
Set<Path> 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<Semver> 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
Expand Down
124 changes: 124 additions & 0 deletions src/java/org/apache/cassandra/utils/FBUtilities.java
Expand Up @@ -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;
Expand All @@ -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;
Expand All @@ -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;
Expand Down Expand Up @@ -131,11 +138,19 @@ public class FBUtilities

private static int availableProcessors = CASSANDRA_AVAILABLE_PROCESSORS.getInt(DatabaseDescriptor.getAvailableProcessors());

private static volatile Supplier<Semver> kernelVersionSupplier = Suppliers.memoize(FBUtilities::getKernelVersionFromUname);

public static void setAvailableProcessors(int value)
{
availableProcessors = value;
}

@VisibleForTesting
public static void setKernelVersionSupplier(Supplier<Semver> supplier)
{
kernelVersionSupplier = supplier;
}

public static int getAvailableProcessors()
{
if (availableProcessors > 0)
Expand Down Expand Up @@ -1080,6 +1095,66 @@ public static void exec(ProcessBuilder pb) throws IOException
}
}

/**
* Starts and waits for the given <code>cmd</code> to finish. If the process does not finish within <code>timeout</code>,
* 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 <code>outBufSize</code>
*/
public static String exec(Map<String, String> 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);
Expand Down Expand Up @@ -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");
}
}

0 comments on commit f0ea12c

Please sign in to comment.