Skip to content

Commit

Permalink
Adds ability to change compaction default in YAML
Browse files Browse the repository at this point in the history
and switches to UCS for the trie unit test suite
  • Loading branch information
blambov committed Apr 20, 2023
1 parent de005f3 commit d27017d
Show file tree
Hide file tree
Showing 5 changed files with 47 additions and 5 deletions.
5 changes: 5 additions & 0 deletions src/java/org/apache/cassandra/config/Config.java
Original file line number Diff line number Diff line change
Expand Up @@ -1087,6 +1087,11 @@ public enum PaxosOnLinearizabilityViolation
public volatile long min_tracked_partition_tombstone_count = 5000;
public volatile boolean top_partitions_enabled = true;

/**
* Default compaction configuration, used if a table does not specify any.
*/
public ParameterizedClass default_compaction = null;

public static Supplier<Config> getOverrideLoadConfig()
{
return overrideLoadConfig;
Expand Down
5 changes: 5 additions & 0 deletions src/java/org/apache/cassandra/config/DatabaseDescriptor.java
Original file line number Diff line number Diff line change
Expand Up @@ -4695,4 +4695,9 @@ public static void setDynamicDataMaskingEnabled(boolean enabled)
conf.dynamic_data_masking_enabled = enabled;
}
}

public static ParameterizedClass getDefaultCompaction()
{
return conf != null ? conf.default_compaction : null;
}
}
21 changes: 19 additions & 2 deletions src/java/org/apache/cassandra/schema/CompactionParams.java
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.config.ParameterizedClass;
import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
import org.apache.cassandra.db.compaction.LeveledCompactionStrategy;
import org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy;
Expand Down Expand Up @@ -84,8 +86,23 @@ public static Optional<TombstoneOption> forName(String name)
ImmutableMap.of(Option.MIN_THRESHOLD.toString(), Integer.toString(DEFAULT_MIN_THRESHOLD),
Option.MAX_THRESHOLD.toString(), Integer.toString(DEFAULT_MAX_THRESHOLD));

public static final CompactionParams DEFAULT =
new CompactionParams(SizeTieredCompactionStrategy.class, DEFAULT_THRESHOLDS, DEFAULT_ENABLED, DEFAULT_PROVIDE_OVERLAPPING_TOMBSTONES);
public static final CompactionParams DEFAULT;
static
{
ParameterizedClass defaultCompaction = DatabaseDescriptor.getDefaultCompaction();
if (defaultCompaction == null)
{
DEFAULT = new CompactionParams(SizeTieredCompactionStrategy.class,
DEFAULT_THRESHOLDS,
DEFAULT_ENABLED,
DEFAULT_PROVIDE_OVERLAPPING_TOMBSTONES);
}
else
{
DEFAULT = create(classFromName(defaultCompaction.class_name),
defaultCompaction.parameters);
}
}

private final Class<? extends AbstractCompactionStrategy> klass;
private final ImmutableMap<String, String> options;
Expand Down
7 changes: 7 additions & 0 deletions test/conf/trie_memtable.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,13 @@
# specific language governing permissions and limitations
# under the License.

# Change default memtable implementation to TrieMemtable
# Note: this attaches at the end of cassandra.yaml, where the memtable configuration setting must be.
default:
inherits: trie

# Change default compaction to UCS
default_compaction:
class_name: UnifiedCompactionStrategy
parameters:
base_shard_count: 1
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.cassandra.cql3.statements;

import java.util.Iterator;
import java.util.Map;
import java.util.Optional;

import com.google.common.collect.ImmutableList;
Expand All @@ -34,9 +35,11 @@

import org.apache.cassandra.config.DatabaseDescriptor;
import org.apache.cassandra.cql3.CQLTester;
import org.apache.cassandra.cql3.CqlBuilder;
import org.apache.cassandra.dht.Token;
import org.apache.cassandra.locator.InetAddressAndPort;
import org.apache.cassandra.locator.TokenMetadata;
import org.apache.cassandra.schema.CompactionParams;
import org.apache.cassandra.schema.Schema;
import org.apache.cassandra.schema.TableId;
import org.apache.cassandra.schema.TableMetadata;
Expand Down Expand Up @@ -896,21 +899,26 @@ private static String tableParametersCql()
" AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'}\n" +
" AND cdc = false\n" +
" AND comment = ''\n" +
" AND compaction = {'class': 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 'max_threshold': '32', 'min_threshold': '4'}\n" +
" AND compaction = " + cqlQuoted(CompactionParams.DEFAULT.asMap()) + "\n" +
" AND compression = {'chunk_length_in_kb': '16', 'class': 'org.apache.cassandra.io.compress.LZ4Compressor'}\n" +
" AND memtable = 'default'\n" +
" AND crc_check_chance = 1.0\n" +
" AND default_time_to_live = 0\n" +
" AND extensions = {}\n" +
" AND gc_grace_seconds = 864000\n" +
" AND incremental_backups = true\n" +
" AND incremental_backups = true\n" +
" AND max_index_interval = 2048\n" +
" AND memtable_flush_period_in_ms = 0\n" +
" AND min_index_interval = 128\n" +
" AND read_repair = 'BLOCKING'\n" +
" AND speculative_retry = '99p';";
}

private static String cqlQuoted(Map<String, String> map)
{
return new CqlBuilder().append(map).toString();
}

private static String mvParametersCql()
{
return "additional_write_policy = '99p'\n" +
Expand All @@ -919,7 +927,7 @@ private static String mvParametersCql()
" AND caching = {'keys': 'ALL', 'rows_per_partition': 'NONE'}\n" +
" AND cdc = false\n" +
" AND comment = ''\n" +
" AND compaction = {'class': 'org.apache.cassandra.db.compaction.SizeTieredCompactionStrategy', 'max_threshold': '32', 'min_threshold': '4'}\n" +
" AND compaction = " + cqlQuoted(CompactionParams.DEFAULT.asMap()) + "\n" +
" AND compression = {'chunk_length_in_kb': '16', 'class': 'org.apache.cassandra.io.compress.LZ4Compressor'}\n" +
" AND memtable = 'default'\n" +
" AND crc_check_chance = 1.0\n" +
Expand Down

0 comments on commit d27017d

Please sign in to comment.