in src/java/org/apache/cassandra/db/guardrails/Guardrails.java [68:557]
isWarning ? format("Creating keyspace %s, current number of keyspaces %s exceeds warning threshold of %s.",
what, value, threshold)
: format("Cannot have more than %s keyspaces, aborting the creation of keyspace %s",
threshold, what));
/**
* Guardrail on the total number of tables on user keyspaces.
*/
public static final MaxThreshold tables =
new MaxThreshold("tables",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getTablesWarnThreshold(),
state -> CONFIG_PROVIDER.getOrCreate(state).getTablesFailThreshold(),
(isWarning, what, value, threshold) ->
isWarning ? format("Creating table %s, current number of tables %s exceeds warning threshold of %s.",
what, value, threshold)
: format("Cannot have more than %s tables, aborting the creation of table %s",
threshold, what));
/**
* Guardrail on the number of columns per table.
*/
public static final MaxThreshold columnsPerTable =
new MaxThreshold("columns_per_table",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getColumnsPerTableWarnThreshold(),
state -> CONFIG_PROVIDER.getOrCreate(state).getColumnsPerTableFailThreshold(),
(isWarning, what, value, threshold) ->
isWarning ? format("The table %s has %s columns, this exceeds the warning threshold of %s.",
what, value, threshold)
: format("Tables cannot have more than %s columns, but %s provided for table %s",
threshold, value, what));
public static final MaxThreshold secondaryIndexesPerTable =
new MaxThreshold("secondary_indexes_per_table",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getSecondaryIndexesPerTableWarnThreshold(),
state -> CONFIG_PROVIDER.getOrCreate(state).getSecondaryIndexesPerTableFailThreshold(),
(isWarning, what, value, threshold) ->
isWarning ? format("Creating secondary index %s, current number of indexes %s exceeds warning threshold of %s.",
what, value, threshold)
: format("Tables cannot have more than %s secondary indexes, aborting the creation of secondary index %s",
threshold, what));
/**
* Guardrail disabling user's ability to create secondary indexes
*/
public static final EnableFlag createSecondaryIndexesEnabled =
new EnableFlag("secondary_indexes",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getSecondaryIndexesEnabled(),
"User creation of secondary indexes");
/**
* Guardrail on the number of materialized views per table.
*/
public static final MaxThreshold materializedViewsPerTable =
new MaxThreshold("materialized_views_per_table",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getMaterializedViewsPerTableWarnThreshold(),
state -> CONFIG_PROVIDER.getOrCreate(state).getMaterializedViewsPerTableFailThreshold(),
(isWarning, what, value, threshold) ->
isWarning ? format("Creating materialized view %s, current number of views %s exceeds warning threshold of %s.",
what, value, threshold)
: format("Tables cannot have more than %s materialized views, aborting the creation of materialized view %s",
threshold, what));
/**
* Guardrail warning about, ignoring or rejecting the usage of certain table properties.
*/
public static final Values<String> tableProperties =
new Values<>("table_properties",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getTablePropertiesWarned(),
state -> CONFIG_PROVIDER.getOrCreate(state).getTablePropertiesIgnored(),
state -> CONFIG_PROVIDER.getOrCreate(state).getTablePropertiesDisallowed(),
"Table Properties");
/**
* Guardrail disabling user-provided timestamps.
*/
public static final EnableFlag userTimestampsEnabled =
new EnableFlag("user_timestamps",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getUserTimestampsEnabled(),
"User provided timestamps (USING TIMESTAMP)");
public static final EnableFlag groupByEnabled =
new EnableFlag("group_by",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getGroupByEnabled(),
"GROUP BY functionality");
/**
* Guardrail disabling ALTER TABLE column mutation access.
*/
public static final EnableFlag alterTableEnabled =
new EnableFlag("alter_table",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getAlterTableEnabled(),
"User access to ALTER TABLE statement for column mutation");
/**
* Guardrail disabling DROP / TRUNCATE TABLE behavior
*/
public static final EnableFlag dropTruncateTableEnabled =
new EnableFlag("drop_truncate_table_enabled",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getDropTruncateTableEnabled(),
"DROP and TRUNCATE TABLE functionality");
/**
* Guardrail disabling DROP KEYSPACE behavior
*/
public static final EnableFlag dropKeyspaceEnabled =
new EnableFlag("drop_keyspace_enabled",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getDropKeyspaceEnabled(),
"DROP KEYSPACE functionality");
/**
* Guardrail disabling bulk loading of SSTables
*/
public static final EnableFlag bulkLoadEnabled =
(EnableFlag) new EnableFlag("bulk_load_enabled",
"Bulk loading of SSTables might potentially destabilize the node.",
state -> CONFIG_PROVIDER.getOrCreate(state).getBulkLoadEnabled(),
"Bulk loading of SSTables").throwOnNullClientState(true);
/**
* Guardrail disabling user's ability to turn off compression
*/
public static final EnableFlag uncompressedTablesEnabled =
new EnableFlag("uncompressed_tables_enabled",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getUncompressedTablesEnabled(),
"Uncompressed table");
/**
* Guardrail disabling the creation of new COMPACT STORAGE tables
*/
public static final EnableFlag compactTablesEnabled =
new EnableFlag("compact_tables",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getCompactTablesEnabled(),
"Creation of new COMPACT STORAGE tables");
/**
* Guardrail to warn or fail a CREATE or ALTER TABLE statement when default_time_to_live is set to 0 and
* the table is using TimeWindowCompactionStrategy compaction or a subclass of it.
*/
public static final EnableFlag zeroTTLOnTWCSEnabled =
new EnableFlag("zero_ttl_on_twcs",
"It is suspicious to use default_time_to_live set to 0 with such compaction strategy. " +
"Please keep in mind that data will not start to automatically expire after they are older " +
"than a respective compaction window unit of a certain size. Please set TTL for your INSERT or UPDATE " +
"statements if you expect data to be expired as table settings will not do it. ",
state -> CONFIG_PROVIDER.getOrCreate(state).getZeroTTLOnTWCSWarned(),
state -> CONFIG_PROVIDER.getOrCreate(state).getZeroTTLOnTWCSEnabled(),
"0 default_time_to_live on a table with " + TimeWindowCompactionStrategy.class.getSimpleName() + " compaction strategy");
/**
* Guardrail to warn on or fail filtering queries that contain intersections on mutable columns at consistency
* levels that require coordinator reconciliation.
*
* @see <a href="https://issues.apache.org/jira/browse/CASSANDRA-19007">CASSANDRA-19007</a>
*/
public static final EnableFlag intersectFilteringQueryEnabled =
new EnableFlag("intersect_filtering_query",
"Filtering queries involving an intersection on multiple mutable (i.e. non-key) columns " +
"over unrepaired data at read consistency levels that would require coordinator " +
"reconciliation may violate the guarantees of those consistency levels.",
state -> CONFIG_PROVIDER.getOrCreate(state).getIntersectFilteringQueryWarned(),
state -> CONFIG_PROVIDER.getOrCreate(state).getIntersectFilteringQueryEnabled(),
"Filtering query with intersection on mutable columns at consistency level requiring coordinator reconciliation");
/**
* Guardrail on the number of elements returned within page.
*/
public static final MaxThreshold pageSize =
new MaxThreshold("page_size",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getPageSizeWarnThreshold(),
state -> CONFIG_PROVIDER.getOrCreate(state).getPageSizeFailThreshold(),
(isWarning, what, value, threshold) ->
isWarning ? format("Query for table %s with page size %s exceeds warning threshold of %s.",
what, value, threshold)
: format("Aborting query for table %s, page size %s exceeds fail threshold of %s.",
what, value, threshold));
/**
* Guardrail on the number of partition keys in the IN clause.
*/
public static final MaxThreshold partitionKeysInSelect =
new MaxThreshold("partition_keys_in_select",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getPartitionKeysInSelectWarnThreshold(),
state -> CONFIG_PROVIDER.getOrCreate(state).getPartitionKeysInSelectFailThreshold(),
(isWarning, what, value, threshold) ->
isWarning ? format("Query with partition keys in IN clause on table %s, with number of " +
"partition keys %s exceeds warning threshold of %s.",
what, value, threshold)
: format("Aborting query with partition keys in IN clause on table %s, " +
"number of partition keys %s exceeds fail threshold of %s.",
what, value, threshold));
/**
* Guardrail disabling operations on lists that require read before write.
*/
public static final EnableFlag readBeforeWriteListOperationsEnabled =
new EnableFlag("read_before_write_list_operations",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getReadBeforeWriteListOperationsEnabled(),
"List operation requiring read before write");
/**
* Guardrail disabling ALLOW FILTERING statement within a query
*/
public static final EnableFlag allowFilteringEnabled =
new EnableFlag("allow_filtering",
"ALLOW FILTERING can potentially visit all the data in the table and have unpredictable performance.",
state -> CONFIG_PROVIDER.getOrCreate(state).getAllowFilteringEnabled(),
"Querying with ALLOW FILTERING");
/**
* Guardrail disabling setting SimpleStrategy via keyspace creation or alteration
*/
public static final EnableFlag simpleStrategyEnabled =
new EnableFlag("simplestrategy",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getSimpleStrategyEnabled(),
"SimpleStrategy");
/**
* Guardrail on the number of restrictions created by a cartesian product of a CQL's {@code IN} query.
*/
public static final MaxThreshold inSelectCartesianProduct =
new MaxThreshold("in_select_cartesian_product",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getInSelectCartesianProductWarnThreshold(),
state -> CONFIG_PROVIDER.getOrCreate(state).getInSelectCartesianProductFailThreshold(),
(isWarning, what, value, threshold) ->
isWarning ? format("The cartesian product of the IN restrictions on %s produces %s values, " +
"this exceeds warning threshold of %s.",
what, value, threshold)
: format("Aborting query because the cartesian product of the IN restrictions on %s " +
"produces %s values, this exceeds fail threshold of %s.",
what, value, threshold));
/**
* Guardrail on read consistency levels.
*/
public static final Values<ConsistencyLevel> readConsistencyLevels =
new Values<>("read_consistency_levels",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getReadConsistencyLevelsWarned(),
state -> Collections.emptySet(),
state -> CONFIG_PROVIDER.getOrCreate(state).getReadConsistencyLevelsDisallowed(),
"read consistency levels");
/**
* Guardrail on write consistency levels.
*/
public static final Values<ConsistencyLevel> writeConsistencyLevels =
new Values<>("write_consistency_levels",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getWriteConsistencyLevelsWarned(),
state -> Collections.emptySet(),
state -> CONFIG_PROVIDER.getOrCreate(state).getWriteConsistencyLevelsDisallowed(),
"write consistency levels");
/**
* Guardrail on the size of a partition.
*/
public static final MaxThreshold partitionSize =
new MaxThreshold("partition_size",
"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 simple type column.
*/
public static final MaxThreshold columnValueSize =
new MaxThreshold("column_value_size",
null,
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getColumnValueSizeWarnThreshold()),
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getColumnValueSizeFailThreshold()),
(isWarning, what, value, threshold) ->
format("Value of column '%s' has size %s, this exceeds the %s threshold of %s.",
what, value, isWarning ? "warning" : "failure", threshold));
/**
* Guardrail on the size of an ascii type column.
*/
public static final MaxThreshold columnAsciiValueSize =
new MaxThreshold("column_ascii_value_size",
null,
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getColumnAsciiValueSizeWarnThreshold()),
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getColumnAsciiValueSizeFailThreshold()),
(isWarning, what, value, threshold) ->
format("Value of column '%s' has size %s, this exceeds the %s threshold of %s.",
what, value, isWarning ? "warning" : "failure", threshold));
/**
* Guardrail on the size of a blob type column.
*/
public static final MaxThreshold columnBlobValueSize =
new MaxThreshold("column_blob_value_size",
null,
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getColumnBlobValueSizeWarnThreshold()),
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getColumnBlobValueSizeFailThreshold()),
(isWarning, what, value, threshold) ->
format("Value of column '%s' has size %s, this exceeds the %s threshold of %s.",
what, value, isWarning ? "warning" : "failure", threshold));
/**
* Guardrail on the size of a text and varchar type column.
*/
public static final MaxThreshold columnTextAndVarcharValueSize =
new MaxThreshold("column_text_value_size",
null,
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getColumnTextAndVarcharValueSizeWarnThreshold()),
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getColumnTextAndVarcharValueSizeFailThreshold()),
(isWarning, what, value, threshold) ->
format("Value of column '%s' has size %s, this exceeds the %s threshold of %s.",
what, value, isWarning ? "warning" : "failure", threshold));
/**
* Guardrail on the size of a collection.
*/
public static final MaxThreshold collectionSize =
new MaxThreshold("collection_size",
null,
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getCollectionSizeWarnThreshold()),
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getCollectionSizeFailThreshold()),
(isWarning, what, value, threshold) ->
format("Detected collection %s of size %s, this exceeds the %s threshold of %s.",
what, value, isWarning ? "warning" : "failure", threshold));
/**
* Guardrail on the size of a map collection.
*/
public static final FallbackThreshold<MaxThreshold> collectionMapSize = new FallbackThreshold<>(
new MaxThreshold("collection_map_size",
null,
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getCollectionMapSizeWarnThreshold()),
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getCollectionMapSizeFailThreshold()),
(isWarning, what, value, threshold) ->
format("Detected collection %s of size %s, this exceeds the %s threshold of %s.",
what, value, isWarning ? "warning" : "failure", threshold)),
collectionSize);
/**
* Guardrail on the size of a set collection.
*/
public static final FallbackThreshold<MaxThreshold> collectionSetSize = new FallbackThreshold<>(
new MaxThreshold("collection_set_size",
null,
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getCollectionSetSizeWarnThreshold()),
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getCollectionSetSizeFailThreshold()),
(isWarning, what, value, threshold) ->
format("Detected collection %s of size %s, this exceeds the %s threshold of %s.",
what, value, isWarning ? "warning" : "failure", threshold))
, collectionSize);
/**
* Guardrail on the size of a list collection.
*/
public static final FallbackThreshold<MaxThreshold> collectionListSize = new FallbackThreshold<>(
new MaxThreshold("collection_list_size",
null,
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getCollectionListSizeWarnThreshold()),
state -> sizeToBytes(CONFIG_PROVIDER.getOrCreate(state).getCollectionListSizeFailThreshold()),
(isWarning, what, value, threshold) ->
format("Detected collection %s of size %s, this exceeds the %s threshold of %s.",
what, value, isWarning ? "warning" : "failure", threshold)),
collectionSize);
/**
* Guardrail on the number of items of a collection.
*/
public static final MaxThreshold itemsPerCollection =
new MaxThreshold("items_per_collection",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getItemsPerCollectionWarnThreshold(),
state -> CONFIG_PROVIDER.getOrCreate(state).getItemsPerCollectionFailThreshold(),
(isWarning, what, value, threshold) ->
format("Detected collection %s with %s items, this exceeds the %s threshold of %s.",
what, value, isWarning ? "warning" : "failure", threshold));
/**
* Guardrail on the number of fields on each UDT.
*/
public static final MaxThreshold fieldsPerUDT =
new MaxThreshold("fields_per_udt",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getFieldsPerUDTWarnThreshold(),
state -> CONFIG_PROVIDER.getOrCreate(state).getFieldsPerUDTFailThreshold(),
(isWarning, what, value, threshold) ->
isWarning ? format("The user type %s has %s columns, this exceeds the warning threshold of %s.",
what, value, threshold)
: format("User types cannot have more than %s columns, but %s provided for user type %s.",
threshold, value, what));
/**
* Guardrail on the usage of vector type.
*
* This may be useful when running clusters that have diverse clients, where some do not support the vector type.
* Clients that do not support vector may fail to establish sessions or fail to read results containing vector
* columns.
*/
public static final EnableFlag vectorTypeEnabled =
new EnableFlag("vector_type_enabled",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getVectorTypeEnabled(),
"usage of the vector type");
/**
* Guardrail on the number of dimensions of vector columns.
*/
public static final MaxThreshold vectorDimensions =
new MaxThreshold("vector_dimensions",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getVectorDimensionsWarnThreshold(),
state -> CONFIG_PROVIDER.getOrCreate(state).getVectorDimensionsFailThreshold(),
(isWarning, what, value, threshold) ->
format("%s has a vector of %s dimensions, this exceeds the %s threshold of %s.",
what, value, isWarning ? "warning" : "failure", threshold));
/**
* Guardrail on the data disk usage on the local node, used by a periodic task to calculate and propagate that status.
* See {@link org.apache.cassandra.service.disk.usage.DiskUsageMonitor} and {@link DiskUsageBroadcaster}.
*/
public static final PercentageThreshold localDataDiskUsage =
new PercentageThreshold("local_data_disk_usage",
null,
state -> CONFIG_PROVIDER.getOrCreate(state).getDataDiskUsagePercentageWarnThreshold(),
state -> CONFIG_PROVIDER.getOrCreate(state).getDataDiskUsagePercentageFailThreshold(),
(isWarning, what, value, threshold) ->
isWarning ? format("Local data disk usage %s(%s) exceeds warning threshold of %s",
value, what, threshold)
: format("Local data disk usage %s(%s) exceeds failure threshold of %s, " +
"will stop accepting writes",
value, what, threshold));
/**
* Guardrail on the data disk usage on replicas, used at write time to verify the status of the involved replicas.
* See {@link org.apache.cassandra.service.disk.usage.DiskUsageMonitor} and {@link DiskUsageBroadcaster}.
*/
public static final Predicates<InetAddressAndPort> replicaDiskUsage =
new Predicates<>("replica_disk_usage",
null,
state -> DiskUsageBroadcaster.instance::isStuffed,
state -> DiskUsageBroadcaster.instance::isFull,
// not using `value` because it represents replica address which should be hidden from client.
(isWarning, value) ->
isWarning ? "Replica disk usage exceeds warning threshold"
: "Write request failed because disk usage exceeds failure threshold");
/**
* Guardrail on passwords for CREATE / ALTER ROLE statements.
*/
public static final PasswordGuardrail password =
new PasswordGuardrail(() -> CONFIG_PROVIDER.getOrCreate(null).getPasswordValidatorConfig());
static
{
// Avoid spamming with notifications about stuffed/full disks
long minNotifyInterval = CassandraRelevantProperties.DISK_USAGE_NOTIFY_INTERVAL_MS.getLong();
localDataDiskUsage.minNotifyIntervalInMs(minNotifyInterval);
replicaDiskUsage.minNotifyIntervalInMs(minNotifyInterval);
}