Skip to content

4.1: CASSANDRA-19703: Ensure prepared_statement INSERT timestamp precedes eviction DELETE #3917

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions CHANGES.txt
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
4.1.9
* Ensure prepared_statement INSERT timestamp precedes eviction DELETE (CASSANDRA-19703)
* Grant permission on keyspaces system_views and system_virtual_schema not possible (CASSANDRA-20171)
* Fix mixed mode paxos ttl commit hang (CASSANDRA-20514)
* Fix paxos mixed mode infinite loop (CASSANDRA-20493)
Expand Down
22 changes: 22 additions & 0 deletions doc/modules/cassandra/pages/cql/cql_singlefile.adoc
Original file line number Diff line number Diff line change
Expand Up @@ -239,6 +239,28 @@ provide values for `LIMIT`, `TIMESTAMP`, and `TTL` clauses. If anonymous
bind markers are used, the names for the query parameters will be
`[limit]`, `[timestamp]`, and `[ttl]`, respectively.

===== Prepared Statement Caching

Prepared Statements are cached by cassandra in-memory using a
https://github.com/ben-manes/caffeine[Caffeine]-managed cache which
can be configured using
xref:managing/configuration/cass_yaml_file.adoc#_prepared_statements_cache_size[`prepared_statements_cache_size`].
The cache is also persisted to the `system.prepared_statements` table
so it can be preloaded into memory on startup.

To ensure optimal performance, it's important to use a bind `<variable>`
for *all non-constant values* in your CQL statements. If you include
literal values directly in the query instead, each variation will be
treated as a unique statement that must be prepared and cached
separately. This will soon overflow the prepared statement cache,
which is small by design.

When the cache reaches its maximum size, older or less frequently
used statements are
https://github.com/ben-manes/caffeine/wiki/Eviction[evicted],
leading to additional overhead as previously prepared statements must
be re-prepared.

[[dataDefinition]]
=== Data Definition

Expand Down
7 changes: 7 additions & 0 deletions src/java/org/apache/cassandra/cql3/QueryHandler.java
Original file line number Diff line number Diff line change
Expand Up @@ -66,6 +66,12 @@ public static class Prepared

public final MD5Digest resultMetadataId;

/**
* Timestamp of when this prepared statement was created. Used in QueryProcessor.preparedStatements cache
* to ensure that the deletion timestamp always succeeds the insert timestamp.
*/
public final long timestamp;

/**
* Contains the CQL statement source if the statement has been "regularly" perpared via
* {@link QueryHandler#prepare(String, ClientState, Map)}.
Expand All @@ -82,6 +88,7 @@ public Prepared(CQLStatement statement, String rawCQLStatement, boolean fullyQua
this.resultMetadataId = ResultSet.ResultMetadata.fromPrepared(statement).getResultMetadataId();
this.fullyQualified = fullyQualified;
this.keyspace = keyspace;
this.timestamp = ClientState.getTimestamp();
}
}
}
72 changes: 55 additions & 17 deletions src/java/org/apache/cassandra/cql3/QueryProcessor.java
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@

import com.github.benmanes.caffeine.cache.Cache;
import com.github.benmanes.caffeine.cache.Caffeine;
import com.github.benmanes.caffeine.cache.RemovalCause;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Predicate;
import com.google.common.collect.*;
Expand Down Expand Up @@ -101,23 +102,22 @@ public class QueryProcessor implements QueryHandler
// counters. Callers of processStatement are responsible for correctly notifying metrics
public static final CQLMetrics metrics = new CQLMetrics();

// Paging size to use when preloading prepared statements.
public static final int PRELOAD_PREPARED_STATEMENTS_FETCH_SIZE = 5000;

// Size of the prepared statement cache in bytes.
public static long PREPARED_STATEMENT_CACHE_SIZE_BYTES = capacityToBytes(DatabaseDescriptor.getPreparedStatementsCacheSizeMiB());

private static final AtomicInteger lastMinuteEvictionsCount = new AtomicInteger(0);

static
{
preparedStatements = Caffeine.newBuilder()
.executor(ImmediateExecutor.INSTANCE)
.maximumWeight(capacityToBytes(DatabaseDescriptor.getPreparedStatementsCacheSizeMiB()))
.maximumWeight(PREPARED_STATEMENT_CACHE_SIZE_BYTES)
.weigher(QueryProcessor::getSizeOfPreparedStatementForCache)
.removalListener((key, prepared, cause) -> {
MD5Digest md5Digest = (MD5Digest) key;
if (cause.wasEvicted())
{
metrics.preparedStatementsEvicted.inc();
lastMinuteEvictionsCount.incrementAndGet();
SystemKeyspace.removePreparedStatement(md5Digest);
}
}).build();
.removalListener((key, prepared, cause) -> evictPreparedStatement(key, cause))
.build();

ScheduledExecutors.scheduledTasks.scheduleAtFixedRate(() -> {
long count = lastMinuteEvictionsCount.getAndSet(0);
Expand All @@ -131,6 +131,16 @@ public class QueryProcessor implements QueryHandler
DatabaseDescriptor.getPreparedStatementsCacheSizeMiB());
}

private static void evictPreparedStatement(MD5Digest key, RemovalCause cause)
{
if (cause.wasEvicted())
{
metrics.preparedStatementsEvicted.inc();
lastMinuteEvictionsCount.incrementAndGet();
SystemKeyspace.removePreparedStatement(key);
}
}

private static long capacityToBytes(long cacheSizeMB)
{
return cacheSizeMB * 1024 * 1024;
Expand All @@ -155,6 +165,12 @@ private enum InternalStateInstance
}

public void preloadPreparedStatements()
{
preloadPreparedStatements(PRELOAD_PREPARED_STATEMENTS_FETCH_SIZE);
}

@VisibleForTesting
public int preloadPreparedStatements(int pageSize)
{
int count = SystemKeyspace.loadPreparedStatements((id, query, keyspace) -> {
try
Expand All @@ -169,17 +185,18 @@ public void preloadPreparedStatements()
// Preload `null` statement for non-fully qualified statements, since it can't be parsed if loaded from cache and will be dropped
if (!prepared.fullyQualified)
preparedStatements.get(computeId(query, null), (ignored_) -> prepared);
return true;
return prepared;
}
catch (RequestValidationException e)
{
JVMStabilityInspector.inspectThrowable(e);
logger.warn(String.format("Prepared statement recreation error, removing statement: %s %s %s", id, query, keyspace));
SystemKeyspace.removePreparedStatement(id);
return false;
return null;
}
});
}, pageSize);
logger.info("Preloaded {} prepared statements", count);
return count;
}


Expand Down Expand Up @@ -556,12 +573,33 @@ public static UntypedResultSet execute(String query, ConsistencyLevel cl, QueryS
public static UntypedResultSet executeInternalWithPaging(String query, int pageSize, Object... values)
{
Prepared prepared = prepareInternal(query);
if (!(prepared.statement instanceof SelectStatement))

return executeInternalWithPaging(prepared.statement, pageSize, values);
}

/**
* Executes with a non-prepared statement using paging. Generally {@link #executeInternalWithPaging(String, int, Object...)}
* should be used instead of this, but this may be used in niche cases like
* {@link SystemKeyspace#loadPreparedStatement(MD5Digest, SystemKeyspace.TriFunction)} where prepared statements are
* being loaded into {@link #preparedStatements} so it doesn't make sense to prepare a statement in this context.
*/
public static UntypedResultSet executeOnceInternalWithPaging(String query, int pageSize, Object... values)
{
QueryState queryState = internalQueryState();
CQLStatement statement = parseStatement(query, queryState.getClientState());
statement.validate(queryState.getClientState());

return executeInternalWithPaging(statement, pageSize, values);
}

private static UntypedResultSet executeInternalWithPaging(CQLStatement statement, int pageSize, Object... values)
{
if (!(statement instanceof SelectStatement))
throw new IllegalArgumentException("Only SELECTs can be paged");

SelectStatement select = (SelectStatement)prepared.statement;
SelectStatement select = (SelectStatement) statement;
int nowInSec = FBUtilities.nowInSeconds();
QueryPager pager = select.getQuery(makeInternalOptionsWithNowInSec(prepared.statement, nowInSec, values), nowInSec).getPager(null, ProtocolVersion.CURRENT);
QueryPager pager = select.getQuery(makeInternalOptionsWithNowInSec(select, nowInSec, values), nowInSec).getPager(null, ProtocolVersion.CURRENT);
return UntypedResultSet.create(select, pager, pageSize);
}

Expand Down Expand Up @@ -800,7 +838,7 @@ public static ResultMessage.Prepared storePreparedStatement(String queryString,

Prepared previous = preparedStatements.get(statementId, (ignored_) -> prepared);
if (previous == prepared)
SystemKeyspace.writePreparedStatement(keyspace, statementId, queryString);
SystemKeyspace.writePreparedStatement(keyspace, statementId, queryString, prepared.timestamp);

ResultSet.PreparedMetadata preparedMetadata = ResultSet.PreparedMetadata.fromPrepared(prepared.statement);
ResultSet.ResultMetadata resultMetadata = ResultSet.ResultMetadata.fromPrepared(prepared.statement);
Expand Down
53 changes: 45 additions & 8 deletions src/java/org/apache/cassandra/db/SystemKeyspace.java
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@
import org.slf4j.LoggerFactory;

import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.cql3.QueryHandler.Prepared;
import org.apache.cassandra.cql3.QueryProcessor;
import org.apache.cassandra.cql3.UntypedResultSet;
import org.apache.cassandra.cql3.functions.AggregateFcts;
Expand Down Expand Up @@ -87,6 +88,7 @@
import org.apache.cassandra.io.util.DataInputBuffer;
import org.apache.cassandra.io.util.DataOutputBuffer;
import org.apache.cassandra.io.util.File;
import org.apache.cassandra.io.util.FileUtils;
import org.apache.cassandra.io.util.RebufferingInputStream;
import org.apache.cassandra.locator.IEndpointSnitch;
import org.apache.cassandra.locator.InetAddressAndPort;
Expand Down Expand Up @@ -130,9 +132,11 @@
import static java.util.concurrent.TimeUnit.MICROSECONDS;
import static org.apache.cassandra.config.Config.PaxosStatePurging.legacy;
import static org.apache.cassandra.config.DatabaseDescriptor.paxosStatePurging;
import static org.apache.cassandra.cql3.QueryProcessor.PREPARED_STATEMENT_CACHE_SIZE_BYTES;
import static org.apache.cassandra.cql3.QueryProcessor.executeInternal;
import static org.apache.cassandra.cql3.QueryProcessor.executeInternalWithNowInSec;
import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternal;
import static org.apache.cassandra.cql3.QueryProcessor.executeOnceInternalWithPaging;
import static org.apache.cassandra.service.paxos.Commit.latest;
import static org.apache.cassandra.utils.CassandraVersion.NULL_VERSION;
import static org.apache.cassandra.utils.CassandraVersion.UNREADABLE_VERSION;
Expand Down Expand Up @@ -1842,11 +1846,11 @@ private static Range<Token> byteBufferToRange(ByteBuffer rawRange, IPartitioner
}
}

public static void writePreparedStatement(String loggedKeyspace, MD5Digest key, String cql)
public static void writePreparedStatement(String loggedKeyspace, MD5Digest key, String cql, long timestamp)
{
executeInternal(format("INSERT INTO %s (logged_keyspace, prepared_id, query_string) VALUES (?, ?, ?)",
executeInternal(format("INSERT INTO %s (logged_keyspace, prepared_id, query_string) VALUES (?, ?, ?) USING TIMESTAMP ?",
PreparedStatements.toString()),
loggedKeyspace, key.byteBuffer(), cql);
loggedKeyspace, key.byteBuffer(), cql, timestamp);
logger.debug("stored prepared statement for logged keyspace '{}': '{}'", loggedKeyspace, cql);
}

Expand All @@ -1862,17 +1866,50 @@ public static void resetPreparedStatements()
preparedStatements.truncateBlockingWithoutSnapshot();
}

public static int loadPreparedStatements(TriFunction<MD5Digest, String, String, Boolean> onLoaded)
public static int loadPreparedStatements(TriFunction<MD5Digest, String, String, Prepared> onLoaded)
{
return loadPreparedStatements(onLoaded, QueryProcessor.PRELOAD_PREPARED_STATEMENTS_FETCH_SIZE);
}

public static int loadPreparedStatements(TriFunction<MD5Digest, String, String, Prepared> onLoaded, int pageSize)
{
String query = String.format("SELECT prepared_id, logged_keyspace, query_string FROM %s.%s", SchemaConstants.SYSTEM_KEYSPACE_NAME, PREPARED_STATEMENTS);
UntypedResultSet resultSet = executeOnceInternal(query);
UntypedResultSet resultSet = executeOnceInternalWithPaging(query, pageSize);
int counter = 0;

// As the cache size may be briefly exceeded before statements are evicted, we allow loading 110% the cache size
// to avoid logging early.
long preparedBytesLoadThreshold = (long) (PREPARED_STATEMENT_CACHE_SIZE_BYTES * 1.1);
long preparedBytesLoaded = 0L;
for (UntypedResultSet.Row row : resultSet)
{
if (onLoaded.accept(MD5Digest.wrap(row.getByteArray("prepared_id")),
row.getString("query_string"),
row.has("logged_keyspace") ? row.getString("logged_keyspace") : null))
Prepared prepared = onLoaded.accept(MD5Digest.wrap(row.getByteArray("prepared_id")),
row.getString("query_string"),
row.has("logged_keyspace") ? row.getString("logged_keyspace") : null);
if (prepared != null)
{
counter++;
preparedBytesLoaded += Math.max(0, prepared.pstmntSize);

if (preparedBytesLoaded > preparedBytesLoadThreshold)
{
// In the event that we detect that we have loaded more bytes than the cache size return early to
// prevent an indefinite startup time. This is almost certainly caused by the prepared statement cache
// leaking (CASSANDRA-19703) which should not recur after being on a version running this code.
// In such a case it's better to warn and continue startup than to continually page over millions of
// prepared statements that would be immediately evicted.
logger.warn("Detected prepared statement cache filling up during preload after preparing {} " +
"statements (loaded {} with prepared_statements_cache_size being {}). " +
"This could be an indication that prepared statements leaked prior to CASSANDRA-19703 " +
"being fixed. Returning early to prevent indefinite startup. " +
"Consider truncating {}.{} to clear out leaked prepared statements.",
counter,
FileUtils.stringifyFileSize(preparedBytesLoaded),
FileUtils.stringifyFileSize(PREPARED_STATEMENT_CACHE_SIZE_BYTES),
SchemaConstants.SYSTEM_KEYSPACE_NAME, PREPARED_STATEMENTS);
break;
}
}
}
return counter;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@
import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
import net.bytebuddy.implementation.MethodDelegation;
import org.apache.cassandra.cql3.CQLStatement;
import org.apache.cassandra.cql3.QueryHandler;
import org.apache.cassandra.cql3.QueryHandler.Prepared;
import org.apache.cassandra.cql3.QueryProcessor;
import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.distributed.api.ConsistencyLevel;
Expand Down Expand Up @@ -268,9 +268,10 @@ public void mixedModeFuzzTest() throws Throwable

c.get(nodeWithFix.get() ? 1 : 2).runOnInstance(() -> {
SystemKeyspace.loadPreparedStatements((id, query, keyspace) -> {
Prepared prepared = QueryProcessor.instance.getPrepared(id);
if (rng.nextBoolean())
QueryProcessor.instance.evictPrepared(id);
return true;
return prepared;
});
});
break;
Expand Down Expand Up @@ -450,7 +451,7 @@ public static ResultMessage.Prepared prepare(String queryString, ClientState cli
if (existing != null)
return existing;

QueryHandler.Prepared prepared = QueryProcessor.parseAndPrepare(queryString, clientState, false);
Prepared prepared = QueryProcessor.parseAndPrepare(queryString, clientState, false);
CQLStatement statement = prepared.statement;

int boundTerms = statement.getBindVariables().size();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
import net.bytebuddy.dynamic.DynamicType;
import net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
import net.bytebuddy.implementation.MethodDelegation;
import org.apache.cassandra.cql3.QueryHandler.Prepared;
import org.apache.cassandra.cql3.QueryProcessor;
import org.apache.cassandra.db.SystemKeyspace;
import org.apache.cassandra.distributed.api.ConsistencyLevel;
Expand Down Expand Up @@ -226,9 +227,10 @@ public void fuzzTest() throws Throwable
case CLEAR_CACHES:
c.get(1).runOnInstance(() -> {
SystemKeyspace.loadPreparedStatements((id, query, keyspace) -> {
Prepared prepared = QueryProcessor.instance.getPrepared(id);
if (rng.nextBoolean())
QueryProcessor.instance.evictPrepared(id);
return true;
return prepared;
});
});
break;
Expand Down
Loading