isWarning ? format()

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);
    }