Skip to content

Commit

Permalink
Add guardrail for partition tombstones and deprecate compaction_tombs…
Browse files Browse the repository at this point in the history
…tone_warning_threshold

patch by Andrés de la Peña; reviewed by Berenguer Blasi and Maxwell Guo for CASSANDRA-17194
  • Loading branch information
adelapena committed Jun 1, 2023
1 parent 4bcadc6 commit b72da02
Show file tree
Hide file tree
Showing 15 changed files with 402 additions and 20 deletions.
1 change: 1 addition & 0 deletions CHANGES.txt
@@ -1,4 +1,5 @@
5.0
* Add guardrail for partition tombstones and deprecate compaction_tombstone_warning_threshold (CASSANDRA-17194)
* Print header and statistics for cassandra-stress output with arbitrary frequency (CASSANDRA-12972)
* CEP-25: Trie-indexed SSTable format (CASSANDRA-18398)
* Make cassandra-stress able to read all credentials from a file (CASSANDRA-18544)
Expand Down
4 changes: 4 additions & 0 deletions NEWS.txt
Expand Up @@ -83,6 +83,7 @@ New features
- Whether DROP KEYSPACE commands are allowed.
- Column value size
- Partition size
- Partition tombstones
- It is possible to list ephemeral snapshots by nodetool listsnaphots command when flag "-e" is specified.
- Added a new flag to `nodetool profileload` and JMX endpoint to set up recurring profile load generation on specified
intervals (see CASSANDRA-17821)
Expand Down Expand Up @@ -193,6 +194,9 @@ Deprecation
- The config property `compaction_large_partition_warning_threshold` has been deprecated in favour of the new
guardrail for partition size. That guardrail is based on the properties `partition_size_warn_threshold` and
`partition_size_fail_threshold`. The warn threshold has a very similar behaviour to the old config property.
- The config property `compaction_tombstone_warning_threshold` has been deprecated in favour of the new guardrail
for partition tombstones. That guardrail is based on the properties `partition_tombstones_warn_threshold` and
`partition_tombstones_fail_threshold`. The warn threshold has a very similar behaviour to the old config property.

4.1
===
Expand Down
12 changes: 11 additions & 1 deletion conf/cassandra.yaml
Expand Up @@ -1564,7 +1564,8 @@ unlogged_batch_across_partitions_warn_threshold: 10
# As of Cassandra 5.0, this property is deprecated in favour of partition_size_warn_threshold.
compaction_large_partition_warning_threshold: 100MiB

# Log a warning when writing more tombstones than this value to a partition
# Log a warning when writing more tombstones than this value to a partition.
# As of Cassandra 5.0, this property is deprecated in favour of partition_tombstones_warn_threshold.
compaction_tombstone_warning_threshold: 100000

# GC Pauses greater than 200 ms will be logged at INFO level
Expand Down Expand Up @@ -1842,6 +1843,15 @@ drop_compact_storage_enabled: false
# partition_size_warn_threshold:
# partition_size_fail_threshold:
#
# Guardrail to warn or fail when writing partitions with more tombstones than threshold.
# The guardrail is only checked when writing sstables (flush and compaction), and exceeding the fail threshold on that
# moment will only log an error message, without interrupting the operation.
# This operates on a per-sstable basis, so it won't detect a large partition if it is spread across multiple sstables.
# The warning threshold replaces the deprecated config property compaction_tombstone_warning_threshold.
# The two thresholds default to -1 to disable.
# partition_tombstones_warn_threshold: -1
# partition_tombstones_fail_threshold: -1
#
# Guardrail to warn or fail when writing column values larger than threshold.
# This guardrail is only applied to the values of regular columns because both the serialized partitions keys and the
# values of the components of the clustering key already have a fixed, relatively small size limit of 65535 bytes, which
Expand Down
3 changes: 3 additions & 0 deletions src/java/org/apache/cassandra/config/Config.java
Expand Up @@ -332,6 +332,7 @@ public MemtableOptions()
public volatile DataStorageSpec.IntMebibytesBound compaction_large_partition_warning_threshold = new DataStorageSpec.IntMebibytesBound("100MiB");
@Replaces(oldName = "min_free_space_per_drive_in_mb", converter = Converters.MEBIBYTES_DATA_STORAGE_INT, deprecated = true)
public DataStorageSpec.IntMebibytesBound min_free_space_per_drive = new DataStorageSpec.IntMebibytesBound("50MiB");
@Deprecated
public volatile Integer compaction_tombstone_warning_threshold = 100000;

// fraction of free disk space available for compaction after min free space is subtracted
Expand Down Expand Up @@ -875,6 +876,8 @@ public static void setClientMode(boolean clientMode)
public volatile boolean simplestrategy_enabled = true;
public volatile DataStorageSpec.LongBytesBound partition_size_warn_threshold = null;
public volatile DataStorageSpec.LongBytesBound partition_size_fail_threshold = null;
public volatile long partition_tombstones_warn_threshold = -1;
public volatile long partition_tombstones_fail_threshold = -1;
public volatile DataStorageSpec.LongBytesBound column_value_size_warn_threshold = null;
public volatile DataStorageSpec.LongBytesBound column_value_size_fail_threshold = null;
public volatile DataStorageSpec.LongBytesBound collection_size_warn_threshold = null;
Expand Down
2 changes: 2 additions & 0 deletions src/java/org/apache/cassandra/config/DatabaseDescriptor.java
Expand Up @@ -2178,11 +2178,13 @@ public static long getCompactionLargePartitionWarningThreshold()
return conf.compaction_large_partition_warning_threshold.toBytesInLong();
}

@Deprecated
public static int getCompactionTombstoneWarningThreshold()
{
return conf.compaction_tombstone_warning_threshold;
}

@Deprecated
public static void setCompactionTombstoneWarningThreshold(int count)
{
conf.compaction_tombstone_warning_threshold = count;
Expand Down
26 changes: 26 additions & 0 deletions src/java/org/apache/cassandra/config/GuardrailsOptions.java
Expand Up @@ -77,6 +77,7 @@ public GuardrailsOptions(Config config)
config.write_consistency_levels_warned = validateConsistencyLevels(config.write_consistency_levels_warned, "write_consistency_levels_warned");
config.write_consistency_levels_disallowed = validateConsistencyLevels(config.write_consistency_levels_disallowed, "write_consistency_levels_disallowed");
validateSizeThreshold(config.partition_size_warn_threshold, config.partition_size_fail_threshold, false, "partition_size");
validateMaxLongThreshold(config.partition_tombstones_warn_threshold, config.partition_tombstones_fail_threshold, "partition_tombstones", false);
validateSizeThreshold(config.column_value_size_warn_threshold, config.column_value_size_fail_threshold, false, "column_value_size");
validateSizeThreshold(config.collection_size_warn_threshold, config.collection_size_fail_threshold, false, "collection_size");
validateMaxIntThreshold(config.items_per_collection_warn_threshold, config.items_per_collection_fail_threshold, "items_per_collection");
Expand Down Expand Up @@ -567,6 +568,31 @@ public void setPartitionSizeThreshold(@Nullable DataStorageSpec.LongBytesBound w
x -> config.partition_size_fail_threshold = x);
}

@Override
public long getPartitionTombstonesWarnThreshold()
{
return config.partition_tombstones_warn_threshold;
}

@Override
public long getPartitionTombstonesFailThreshold()
{
return config.partition_tombstones_fail_threshold;
}

public void setPartitionTombstonesThreshold(long warn, long fail)
{
validateMaxLongThreshold(warn, fail, "partition_tombstones", false);
updatePropertyWithLogging("partition_tombstones_warn_threshold",
warn,
() -> config.partition_tombstones_warn_threshold,
x -> config.partition_tombstones_warn_threshold = x);
updatePropertyWithLogging("partition_tombstones_fail_threshold",
fail,
() -> config.partition_tombstones_fail_threshold,
x -> config.partition_tombstones_fail_threshold = x);
}

@Override
@Nullable
public DataStorageSpec.LongBytesBound getColumnValueSizeWarnThreshold()
Expand Down
32 changes: 31 additions & 1 deletion src/java/org/apache/cassandra/db/guardrails/Guardrails.java
Expand Up @@ -316,13 +316,25 @@ public final class Guardrails implements GuardrailsMBean
*/
public static final MaxThreshold partitionSize =
new MaxThreshold("partition_size",
"Too large partitions can cause performance problems. ",
"Too large partitions can cause performance problems.",
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getPartitionSizeWarnThreshold()),
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getPartitionSizeFailThreshold()),
(isWarning, what, value, threshold) ->
format("Partition %s has size %s, this exceeds the %s threshold of %s.",
what, value, isWarning ? "warning" : "failure", threshold));

/**
* Guardrail on the number of rows of a partition.
*/
public static final MaxThreshold partitionTombstones =
new MaxThreshold("partition_tombstones",
"Partitions with too many tombstones can cause performance problems.",
state -> CONFIG_PROVIDER.getOrCreate(state).getPartitionTombstonesWarnThreshold(),
state -> CONFIG_PROVIDER.getOrCreate(state).getPartitionTombstonesFailThreshold(),
(isWarning, what, value, threshold) ->
format("Partition %s has %s tombstones, this exceeds the %s threshold of %s.",
what, value, isWarning ? "warning" : "failure", threshold));

/**
* Guardrail on the size of a collection.
*/
Expand Down Expand Up @@ -823,6 +835,24 @@ public void setPartitionSizeThreshold(@Nullable String warnSize, @Nullable Strin
DEFAULT_CONFIG.setPartitionSizeThreshold(sizeFromString(warnSize), sizeFromString(failSize));
}

@Override
public long getPartitionTombstonesWarnThreshold()
{
return DEFAULT_CONFIG.getPartitionTombstonesWarnThreshold();
}

@Override
public long getPartitionTombstonesFailThreshold()
{
return DEFAULT_CONFIG.getPartitionTombstonesFailThreshold();
}

@Override
public void setPartitionTombstonesThreshold(long warn, long fail)
{
DEFAULT_CONFIG.setPartitionTombstonesThreshold(warn, fail);
}

@Override
@Nullable
public String getColumnValueSizeWarnThreshold()
Expand Down
10 changes: 10 additions & 0 deletions src/java/org/apache/cassandra/db/guardrails/GuardrailsConfig.java
Expand Up @@ -250,6 +250,16 @@ public interface GuardrailsConfig
@Nullable
DataStorageSpec.LongBytesBound getPartitionSizeFailThreshold();

/**
* @return The threshold to warn when writing partitions with more tombstones than threshold.
*/
long getPartitionTombstonesWarnThreshold();

/**
* @return The threshold to fail when writing partitions with more tombstones than threshold.
*/
long getPartitionTombstonesFailThreshold();

/**
* @return The threshold to warn when writing column values larger than threshold.
*/
Expand Down
26 changes: 23 additions & 3 deletions src/java/org/apache/cassandra/db/guardrails/GuardrailsMBean.java
Expand Up @@ -372,21 +372,21 @@ public interface GuardrailsMBean
* @return The threshold to warn when an IN query creates a cartesian product with a size exceeding threshold.
* -1 means disabled.
*/
public int getInSelectCartesianProductWarnThreshold();
int getInSelectCartesianProductWarnThreshold();

/**
* @return The threshold to prevent IN queries creating a cartesian product with a size exceeding threshold.
* -1 means disabled.
*/
public int getInSelectCartesianProductFailThreshold();
int getInSelectCartesianProductFailThreshold();

/**
* @param warn The threshold to warn when an IN query creates a cartesian product with a size exceeding threshold.
* -1 means disabled.
* @param fail The threshold to prevent IN queries creating a cartesian product with a size exceeding threshold.
* -1 means disabled.
*/
public void setInSelectCartesianProductThreshold(int warn, int fail);
void setInSelectCartesianProductThreshold(int warn, int fail);

/**
* @return consistency levels that are warned about when reading.
Expand Down Expand Up @@ -495,6 +495,26 @@ public interface GuardrailsMBean
*/
void setPartitionSizeThreshold(@Nullable String warnSize, @Nullable String failSize);

/**
* @return The threshold to warn when encountering partitions with more tombstones than threshold. -1 means disabled.
*/
long getPartitionTombstonesWarnThreshold();

/**
* @return The threshold to fail when encountering partitions with more tombstones than threshold. -1 means disabled.
* Triggering a failure emits a log message and a diagnostic event, but it doesn't throw an exception interrupting
* the offending sstable write.
*/
long getPartitionTombstonesFailThreshold();

/**
* @param warn The threshold to warn when encountering partitions with more tombstones than threshold. -1 means disabled.
* @param fail The threshold to fail when encountering partitions with more tombstones than threshold. -1 means disabled.
* Triggering a failure emits a log message and a diagnostic event, but it desn't throw an exception
* interrupting the offending sstable write.
*/
void setPartitionTombstonesThreshold(long warn, long fail);

/**
* @return The threshold to warn when encountering column values larger than threshold, as a string formatted as
* in, for example, {@code 10GiB}, {@code 20MiB}, {@code 30KiB} or {@code 40B}. A {@code null} value means disabled.
Expand Down
Expand Up @@ -32,6 +32,7 @@
import org.apache.cassandra.db.DeletionPurger;
import org.apache.cassandra.db.DeletionTime;
import org.apache.cassandra.db.guardrails.Guardrails;
import org.apache.cassandra.db.guardrails.Threshold;
import org.apache.cassandra.db.lifecycle.LifecycleNewTracker;
import org.apache.cassandra.db.rows.ComplexColumnData;
import org.apache.cassandra.db.rows.PartitionSerializationException;
Expand Down Expand Up @@ -213,7 +214,8 @@ private AbstractRowIndexEntry endPartition(DecoratedKey key, DeletionTime partit

long endPosition = dataWriter.position();
long rowSize = endPosition - partitionWriter.getInitialPosition();
guardPartitionSize(key, rowSize);
guardPartitionThreshold(Guardrails.partitionSize, key, rowSize);
guardPartitionThreshold(Guardrails.partitionTombstones, key, metadataCollector.totalTombstones);
maybeLogLargePartitionWarning(key, rowSize);
maybeLogManyTombstonesWarning(key, metadataCollector.totalTombstones);
metadataCollector.addPartitionSizeInBytes(rowSize);
Expand Down Expand Up @@ -325,19 +327,20 @@ protected FileHandle openDataFile(long lengthOverride, StatsMetadata statsMetada
return dataFile;
}

private void guardPartitionSize(DecoratedKey key, long rowSize)
private void guardPartitionThreshold(Threshold guardrail, DecoratedKey key, long size)
{
if (Guardrails.partitionSize.triggersOn(rowSize, null))
if (guardrail.triggersOn(size, null))
{
String what = String.format("%s.%s:%s on sstable %s",
metadata.keyspace,
metadata.name,
metadata().partitionKeyType.getString(key.getKey()),
getFilename());
Guardrails.partitionSize.guard(rowSize, what, true, null);
String message = String.format("%s.%s:%s on sstable %s",
metadata.keyspace,
metadata.name,
metadata().partitionKeyType.getString(key.getKey()),
getFilename());
guardrail.guard(size, message, true, null);
}
}

@Deprecated
private void maybeLogLargePartitionWarning(DecoratedKey key, long rowSize)
{
if (rowSize > DatabaseDescriptor.getCompactionLargePartitionWarningThreshold())
Expand All @@ -347,6 +350,7 @@ private void maybeLogLargePartitionWarning(DecoratedKey key, long rowSize)
}
}

@Deprecated
private void maybeLogManyTombstonesWarning(DecoratedKey key, int tombstoneCount)
{
if (tombstoneCount > DatabaseDescriptor.getCompactionTombstoneWarningThreshold())
Expand Down
2 changes: 2 additions & 0 deletions src/java/org/apache/cassandra/service/StorageService.java
Expand Up @@ -6949,6 +6949,7 @@ public void setKeyspaceCountWarnThreshold(int value)
DatabaseDescriptor.setKeyspaceCountWarnThreshold(value);
}

@Override
public void setCompactionTombstoneWarningThreshold(int count)
{
if (count < 0)
Expand All @@ -6957,6 +6958,7 @@ public void setCompactionTombstoneWarningThreshold(int count)
DatabaseDescriptor.setCompactionTombstoneWarningThreshold(count);
}

@Override
public int getCompactionTombstoneWarningThreshold()
{
return DatabaseDescriptor.getCompactionTombstoneWarningThreshold();
Expand Down
Expand Up @@ -1115,8 +1115,10 @@ public void enableAuditLog(String loggerName, String includedKeyspaces, String e
@Deprecated
void setKeyspaceCountWarnThreshold(int value);

public void setCompactionTombstoneWarningThreshold(int count);
public int getCompactionTombstoneWarningThreshold();
@Deprecated
void setCompactionTombstoneWarningThreshold(int count);
@Deprecated
int getCompactionTombstoneWarningThreshold();

public boolean getReadThresholdsEnabled();
public void setReadThresholdsEnabled(boolean value);
Expand Down
Expand Up @@ -75,12 +75,11 @@ protected Cluster getCluster()
@Test
public void testPartitionSize()
{
// test yaml-loaded config
testPartitionSize(WARN_THRESHOLD, FAIL_THRESHOLD);
}
schemaChange("DROP TABLE %s");

@Test
public void testPartitionSizeWithDynamicUpdate()
{
// test dynamic config
int warn = WARN_THRESHOLD * 2;
int fail = FAIL_THRESHOLD * 2;
cluster.get(1).runOnInstance(() -> Guardrails.instance.setPartitionSizeThreshold(warn + "B", fail + "B"));
Expand Down

0 comments on commit b72da02

Please sign in to comment.